eacf7b
@@ -19,6 +19,9 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 import java.io.IOException;
 import java.util.AbstractMap.SimpleEntry;
 import java.util.ArrayList;
@@ -50,8 +53,6 @@
import org.apache.hadoop.hbase.client.AsyncProcess.AsyncRequestFuture;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
 /**
  * HTableMultiplexer provides a thread-safe non blocking PUT API across all the tables.
  * Each put will be sharded into different buffer queues based on its destination region server.
@@ -97,7 +98,18 @@
public class HTableMultiplexer {
    */
   public HTableMultiplexer(Configuration conf, int perRegionServerBufferQueueSize)
       throws IOException {
-    this.conn = (ClusterConnection) ConnectionFactory.createConnection(conf);
+    this(ConnectionFactory.createConnection(conf), conf, perRegionServerBufferQueueSize);
+  }
+
+  /**
+   * @param conn The HBase connection.
+   * @param conf The HBase configuration
+   * @param perRegionServerBufferQueueSize determines the max number of the buffered Put ops for
+   *          each region server before dropping the request.
+   */
+  public HTableMultiplexer(Connection conn, Configuration conf,
+      int perRegionServerBufferQueueSize) {
+    this.conn = (ClusterConnection) conn;
     this.pool = HTable.getDefaultExecutor(conf);
     // how many times we could try in total, one more than retry number
     this.maxAttempts = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
@@ -116,6 +128,18 @@
public class HTableMultiplexer {
     this.workerConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0);
   }
 
+  /**
+   * Closes the internal {@link Connection}. Does nothing if the {@link Connection} has already
+   * been closed.
+   * @throws IOException If there is an error closing the connection.
+   */
+  @SuppressWarnings("deprecation")
+  public synchronized void close() throws IOException {
+    if (!getConnection().isClosed()) {
+      getConnection().close();
+    }
+  }
+
   /**
    * The put request will be buffered by its corresponding buffer queue. Return false if the queue
    * is already full.
@@ -170,13 +194,28 @@
public class HTableMultiplexer {
    * @return true if the request can be accepted by its corresponding buffer queue.
    */
   public boolean put(final TableName tableName, final Put put, int maxAttempts) {
+    return _put(tableName, put, maxAttempts, false);
+  }
+
+  /**
+   * Internal "put" which exposes a boolean flag to control whether or not the region location
+   * cache should be reloaded when trying to queue the {@link Put}.
+   * @param tableName Destination table for the Put
+   * @param put The Put to send
+   * @param maxAttempts Number of attempts to retry the {@code put}
+   * @param reloadCache Should the region location cache be reloaded
+   * @return true if the request was accepted in the queue, otherwise false
+   */
+  boolean _put(final TableName tableName, final Put put, int maxAttempts, boolean reloadCache) {
     if (maxAttempts <= 0) {
       return false;
     }
 
     try {
       HTable.validatePut(put, maxKeyValueSize);
-      HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), false);
+      // Allow mocking to get at the connection, but don't expose the connection to users.
+      ClusterConnection conn = (ClusterConnection) getConnection();
+      HRegionLocation loc = conn.getRegionLocation(tableName, put.getRow(), reloadCache);
       if (loc != null) {
         // Add the put pair into its corresponding queue.
         LinkedBlockingQueue<PutStatus> queue = getQueue(loc);
@@ -215,7 +254,8 @@
public class HTableMultiplexer {
     return new HTableMultiplexerStatus(serverToFlushWorkerMap);
   }
 
-  private LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
+  @VisibleForTesting
+  LinkedBlockingQueue<PutStatus> getQueue(HRegionLocation addr) {
     FlushWorker worker = serverToFlushWorkerMap.get(addr);
     if (worker == null) {
       synchronized (this.serverToFlushWorkerMap) {
@@ -232,6 +272,11 @@
public class HTableMultiplexer {
     return worker.getQueue();
   }
 
+  @VisibleForTesting
+  ClusterConnection getConnection() {
+    return this.conn;
+  }
+
   /**
    * HTableMultiplexerStatus keeps track of the current status of the HTableMultiplexer.
    * report the number of buffered requests and the number of the failed (dropped) requests
@@ -340,10 +385,11 @@
public class HTableMultiplexer {
     }
   }
 
-  private static class PutStatus {
-    private final HRegionInfo regionInfo;
-    private final Put put;
-    private final int maxAttempCount;
+  @VisibleForTesting
+  static class PutStatus {
+    final HRegionInfo regionInfo;
+    final Put put;
+    final int maxAttempCount;
 
     public PutStatus(HRegionInfo regionInfo, Put put, int maxAttempCount) {
       this.regionInfo = regionInfo;
@@ -392,7 +438,8 @@
public class HTableMultiplexer {
     }
   }
 
-  private static class FlushWorker implements Runnable {
+  @VisibleForTesting
+  static class FlushWorker implements Runnable {
     private final HRegionLocation addr;
     private final LinkedBlockingQueue<PutStatus> queue;
     private final HTableMultiplexer multiplexer;
@@ -440,7 +487,7 @@
public class HTableMultiplexer {
       return this.maxLatency.getAndSet(0);
     }
 
-    private boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException {
+    boolean resubmitFailedPut(PutStatus ps, HRegionLocation oldLoc) throws IOException {
       // Decrease the retry count
       final int retryCount = ps.maxAttempCount - 1;
 
@@ -449,10 +496,10 @@
public class HTableMultiplexer {
         return false;
       }
 
-      int cnt = retryInQueue.incrementAndGet();
-      if (cnt > maxRetryInQueue) {
+      int cnt = getRetryInQueue().incrementAndGet();
+      if (cnt > getMaxRetryInQueue()) {
         // Too many Puts in queue for resubmit, give up this
-        retryInQueue.decrementAndGet();
+        getRetryInQueue().decrementAndGet();
         return false;
       }
 
@@ -460,22 +507,21 @@
public class HTableMultiplexer {
       // The currentPut is failed. So get the table name for the currentPut.
       final TableName tableName = ps.regionInfo.getTable();
 
-      long delayMs = ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
-        multiplexer.maxAttempts - retryCount - 1);
+      long delayMs = getNextDelay(retryCount);
       if (LOG.isDebugEnabled()) {
         LOG.debug("resubmitting after " + delayMs + "ms: " + retryCount);
       }
 
-      executor.schedule(new Runnable() {
+      getExecutor().schedule(new Runnable() {
         @Override
         public void run() {
           boolean succ = false;
           try {
-            succ = FlushWorker.this.multiplexer.put(tableName, failedPut, retryCount);
+            succ = FlushWorker.this.getMultiplexer()._put(tableName, failedPut, retryCount, true);
           } finally {
-            FlushWorker.this.retryInQueue.decrementAndGet();
+            FlushWorker.this.getRetryInQueue().decrementAndGet();
             if (!succ) {
-              FlushWorker.this.totalFailedPutCount.incrementAndGet();
+              FlushWorker.this.getTotalFailedPutCount().incrementAndGet();
             }
           }
         }
@@ -483,6 +529,37 @@
public class HTableMultiplexer {
       return true;
     }
 
+    @VisibleForTesting
+    long getNextDelay(int retryCount) {
+      return ConnectionUtils.getPauseTime(multiplexer.flushPeriod,
+          multiplexer.maxAttempts - retryCount - 1);
+    }
+
+    @VisibleForTesting
+    AtomicInteger getRetryInQueue() {
+      return this.retryInQueue;
+    }
+
+    @VisibleForTesting
+    int getMaxRetryInQueue() {
+      return this.maxRetryInQueue;
+    }
+
+    @VisibleForTesting
+    AtomicLong getTotalFailedPutCount() {
+      return this.totalFailedPutCount;
+    }
+
+    @VisibleForTesting
+    HTableMultiplexer getMultiplexer() {
+      return this.multiplexer;
+    }
+
+    @VisibleForTesting
+    ScheduledExecutorService getExecutor() {
+      return this.executor;
+    }
+
     @Override
     public void run() {
       int failedCount = 0;
